[SPARK-16814][SQL] Fix deprecated parquet constructor usage#14419
[SPARK-16814][SQL] Fix deprecated parquet constructor usage#14419holdenk wants to merge 11 commits intoapache:masterfrom
Conversation
|
Test build #63039 has finished for PR 14419 at commit
|
|
Do we know if this works across versions of Parquet that are still in play for Spark builds? I know we've got some YARN and Scala warnings that we can't adjust for that reason. Just double checking. @HyukjinKwon did you look at this too or am I imagining it? |
|
Thank you for cc me! Yes, actually I took a look for this as well and it seems okay (I was thinking doing this with other issues together). |
|
Could I cc @liancheng who has more insight about this please? |
|
So yes as @HyukjinKwon says its just in the tests. |
|
@HyukjinKwon do you have a PR for this? If so I can go ahead and close this one (and lets maybe sync some on the deprecation stuff we are both cleaning up :)) |
|
No, I don't :). I just meant I wanted to say that this fix is reasonable and the reason I didn't submit a PR has not been because there is potential incompatability. It's just been because just I was thinking doing this with other ones in the future. I like this PR :). Sorry for confusing you. |
| val messageType = MessageTypeParser.parseMessageType(schema) | ||
| val writeSupport = new DirectWriteSupport(messageType, metadata) | ||
| val parquetWriter = new ParquetWriter[RecordConsumer => Unit](new Path(path), writeSupport) | ||
| val testWriteSupport = new DirectWriteSupport(messageType, metadata) |
There was a problem hiding this comment.
can you explain what's going on here? This seems like pretty complicated.
There was a problem hiding this comment.
Sure, so the parquetWriter constructors are deprecated now and its been replaced with a builder interface. For Avro and others there is a standard builder - but for sort "raw" formats you need to implement your own builder. This is equivalent to the old constructor we were using - you can see the deprecation in https://github.com/apache/parquet-mr/pull/199/files as well as how the builder interface ends up calling an equivalent (now protected) constructor. Also since our WriteSupport doesn't need to change based on the configuration we always return the same writesupport regardless of conf.
If it would be useful I can add some of this as a comment in the sourcecode.
There was a problem hiding this comment.
@rxin Does this explanation make sense for you?
There was a problem hiding this comment.
You might just go ahead and add it as a comment for good measure.
Isn't @Override the Java annotation? thought Scala needed @override but could be missing something.
There was a problem hiding this comment.
I didn't know Java annotation passes Scala style checking. Should we consider adding a rule for this?
There was a problem hiding this comment.
Sure, because I don't think it would actually cause the Scala compiler to verify it overrides. It's not illegal to use a Java annotation, just doesn't do anything in this case?
There was a problem hiding this comment.
Filed here, https://issues.apache.org/jira/browse/SPARK-16877 but I will do some researches and take a look into this deeper first before opening a PR.
There was a problem hiding this comment.
Ack yeah that's what I meant, thank you.
…es since we just expose getWriteSupport)
|
Test build #63051 has finished for PR 14419 at commit
|
|
@HyukjinKwon no worries - just wanted to make sure we weren't accidentally duplicating eachothers efforts :) |
| val writeSupport = new DirectWriteSupport(messageType, metadata) | ||
| val parquetWriter = new ParquetWriter[RecordConsumer => Unit](new Path(path), writeSupport) | ||
| val testWriteSupport = new DirectWriteSupport(messageType, metadata) | ||
| case class ParquetWriterBuilder() extends |
There was a problem hiding this comment.
One thing I am a bit wondering though is why this has to be case class instead of just class. Because it seems it is self-contained, so it seems not have to be serializable and it seems there is no pattern matching with this. Just curious.
There was a problem hiding this comment.
Thats a good point, they probably don't need to be case classes (just happened to write that way when I started the code) - I'll switch them to regular classes.
…f github.com:holdenk/spark into SPARK-16814-fix-deprecated-parquet-constructor-usage
|
Test build #63062 has finished for PR 14419 at commit
|
| val testWriteSupport = new DirectWriteSupport(messageType, metadata) | ||
| class ParquetWriterBuilder() extends | ||
| ParquetWriter.Builder[RecordConsumer => Unit, ParquetWriterBuilder](new Path(path)) { | ||
| @Override def getWriteSupport(conf: org.apache.hadoop.conf.Configuration) = { |
There was a problem hiding this comment.
Import Configuration? and maybe just write the trivial body of both methods inline without braces on one line. Just personal taste really, to keep this anonymous class compact
|
Test build #63183 has finished for PR 14419 at commit
|
|
Merged to master |
What changes were proposed in this pull request?
Replace deprecated ParquetWriter with the new builders
How was this patch tested?
Existing tests